1   /*
2    * Licensed to the Apache Software Foundation (ASF) under one or more
3    * contributor license agreements.  See the NOTICE file distributed with
4    * this work for additional information regarding copyright ownership.
5    * The ASF licenses this file to You under the Apache License, Version 2.0
6    * (the "License"); you may not use this file except in compliance with
7    * the License.  You may obtain a copy of the License at
8    *
9    *     http://www.apache.org/licenses/LICENSE-2.0
10   *
11   * Unless required by applicable law or agreed to in writing, software
12   * distributed under the License is distributed on an "AS IS" BASIS,
13   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14   * See the License for the specific language governing permissions and
15   * limitations under the License.
16   */
17  
18  package org.apache.solr.search;
19  
20  import java.io.IOException;
21  import java.util.ArrayList;
22  import java.util.Arrays;
23  import java.util.EnumSet;
24  import java.util.HashMap;
25  import java.util.Iterator;
26  import java.util.List;
27  import java.util.Map;
28  
29  import com.carrotsearch.hppc.FloatArrayList;
30  import com.carrotsearch.hppc.IntArrayList;
31  import com.carrotsearch.hppc.IntIntHashMap;
32  import com.carrotsearch.hppc.IntLongHashMap;
33  import com.carrotsearch.hppc.cursors.IntIntCursor;
34  import com.carrotsearch.hppc.cursors.IntLongCursor;
35  
36  import org.apache.commons.lang.StringUtils;
37  
38  import org.apache.lucene.index.DocValues;
39  import org.apache.lucene.index.DocValuesType;
40  import org.apache.lucene.index.FieldInfo;
41  import org.apache.lucene.index.FieldInfos;
42  import org.apache.lucene.index.FilterLeafReader;
43  import org.apache.lucene.index.LeafReader;
44  import org.apache.lucene.index.LeafReaderContext;
45  import org.apache.lucene.index.MultiDocValues;
46  import org.apache.lucene.index.NumericDocValues;
47  import org.apache.lucene.index.SortedDocValues;
48  import org.apache.lucene.queries.function.FunctionQuery;
49  import org.apache.lucene.queries.function.FunctionValues;
50  import org.apache.lucene.queries.function.ValueSource;
51  import org.apache.lucene.search.DocIdSetIterator;
52  import org.apache.lucene.search.FieldComparator;
53  import org.apache.lucene.search.LeafFieldComparator;
54  import org.apache.lucene.search.IndexSearcher;
55  import org.apache.lucene.search.Query;
56  import org.apache.lucene.search.Scorer;
57  import org.apache.lucene.search.Sort;
58  import org.apache.lucene.search.SortField;
59  import org.apache.lucene.uninverting.UninvertingReader;
60  import org.apache.lucene.util.ArrayUtil;
61  import org.apache.lucene.util.BitSetIterator;
62  import org.apache.lucene.util.BytesRef;
63  import org.apache.lucene.util.FixedBitSet;
64  import org.apache.lucene.util.LongValues;
65  import org.apache.solr.common.SolrException;
66  import org.apache.solr.common.params.ModifiableSolrParams;
67  import org.apache.solr.common.params.SolrParams;
68  import org.apache.solr.common.util.NamedList;
69  import org.apache.solr.handler.component.ResponseBuilder;
70  import org.apache.solr.handler.component.QueryElevationComponent;
71  import org.apache.solr.request.LocalSolrQueryRequest;
72  import org.apache.solr.request.SolrQueryRequest;
73  import org.apache.solr.request.SolrRequestInfo;
74  import org.apache.solr.schema.FieldType;
75  import org.apache.solr.schema.StrField;
76  import org.apache.solr.schema.TrieFloatField;
77  import org.apache.solr.schema.TrieIntField;
78  import org.apache.solr.schema.TrieLongField;
79  
80  /**
81  
82   The <b>CollapsingQParserPlugin</b> is a PostFilter that performs field collapsing.
83   This is a high performance alternative to standard Solr
84   field collapsing (with ngroups) when the number of distinct groups
85   in the result set is high.
86   <p>
87   Sample syntax:
88   <p>
89   Collapse based on the highest scoring document:
90   <p>
91  
92   fq=(!collapse field=field_name}
93  
94   <p>
95   Collapse based on the min value of a numeric field:
96   <p>
97   fq={!collapse field=field_name min=field_name}
98   <p>
99   Collapse based on the max value of a numeric field:
100  <p>
101  fq={!collapse field=field_name max=field_name}
102  <p>
103  Collapse with a null policy:
104  <p>
105  fq={!collapse field=field_name nullPolicy=nullPolicy}
106  <p>
107  There are three null policies: <br>
108  ignore : removes docs with a null value in the collapse field (default).<br>
109  expand : treats each doc with a null value in the collapse field as a separate group.<br>
110  collapse : collapses all docs with a null value into a single group using either highest score, or min/max.
111  <p>
112  The CollapsingQParserPlugin fully supports the QueryElevationComponent
113  **/
114 
115 public class CollapsingQParserPlugin extends QParserPlugin {
116 
117   public static final String NAME = "collapse";
118   public static final String NULL_COLLAPSE = "collapse";
119   public static final String NULL_IGNORE = "ignore";
120   public static final String NULL_EXPAND = "expand";
121   public static final String HINT_TOP_FC = "top_fc";
122   public static final String HINT_MULTI_DOCVALUES = "multi_docvalues";
123 
124 
125   public void init(NamedList namedList) {
126 
127   }
128 
129   public QParser createParser(String qstr, SolrParams localParams, SolrParams params, SolrQueryRequest request) {
130     return new CollapsingQParser(qstr, localParams, params, request);
131   }
132 
133   private static class CollapsingQParser extends QParser {
134 
135     public CollapsingQParser(String qstr, SolrParams localParams, SolrParams params, SolrQueryRequest request) {
136       super(qstr, localParams, params, request);
137     }
138 
139     public Query parse() throws SyntaxError {
140       try {
141         return new CollapsingPostFilter(localParams, params, req);
142       } catch (Exception e) {
143         throw new SyntaxError(e.getMessage(), e);
144       }
145     }
146   }
147 
148   public static enum GroupHeadSelectorType {
149     MIN, MAX, SORT, SCORE;
150     public static EnumSet<GroupHeadSelectorType> MIN_MAX = EnumSet.of(MIN, MAX);
151   }
152   
153   /**
154    * Models all the information about how group head documents should be selected
155    */
156   public static final class GroupHeadSelector {
157     
158     /** 
159      * The param value for this selector whose meaning depends on type.  
160      * (ie: a field or valuesource for MIN/MAX, a sort string for SORT, "score" for SCORE). 
161      * Will never be null.
162      */
163     public final String selectorText;
164     /** The type for this selector, will never be null */
165     public final GroupHeadSelectorType type;
166     private GroupHeadSelector(String s, GroupHeadSelectorType type) {
167       assert null != s;
168       assert null != type;
169       
170       this.selectorText = s;
171       this.type = type;
172     }
173 
174     @Override
175     public boolean equals(final Object other) {
176       if (other instanceof GroupHeadSelector) {
177         final GroupHeadSelector that = (GroupHeadSelector) other;
178         return (this.type == that.type) && this.selectorText.equals(that.selectorText);
179       }
180       return false;
181     }
182     
183     @Override
184     public int hashCode() {
185       return 17 * (31 + selectorText.hashCode()) * (31 + type.hashCode());
186     }
187     
188     /**
189      * returns a new GroupHeadSelector based on the specified local params
190      */
191     public static GroupHeadSelector build(final SolrParams localParams) {
192       final String sortString = StringUtils.defaultIfBlank(localParams.get("sort"), null);
193       final String max = StringUtils.defaultIfBlank(localParams.get("max"), null);
194       final String min = StringUtils.defaultIfBlank(localParams.get("min"), null);
195 
196       if (1 < numNotNull(min, max, sortString)) {
197         throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
198                                 "At most one localparam for selecting documents (min, max, sort) may be specified: " + localParams.toString());
199       }
200       
201       if (null != sortString) {
202         return new GroupHeadSelector(sortString, GroupHeadSelectorType.SORT);
203       } else if (null != min) {
204         return new GroupHeadSelector(min, GroupHeadSelectorType.MIN);
205       } else if (null != max) {
206         return new GroupHeadSelector(max, GroupHeadSelectorType.MAX);
207       }
208       // default
209       return new GroupHeadSelector("score", GroupHeadSelectorType.SCORE);
210     }
211   }
212 
213   public static class CollapsingPostFilter extends ExtendedQueryBase implements PostFilter, ScoreFilter {
214 
215     private String collapseField;
216     private final GroupHeadSelector groupHeadSelector;
217     private final SortSpec sortSpec; // may be null, parsed at most once from groupHeadSelector
218     public String hint;
219     private boolean needsScores = true;
220     private int nullPolicy;
221     private Map<BytesRef, Integer> boosted;
222     public static final int NULL_POLICY_IGNORE = 0;
223     public static final int NULL_POLICY_COLLAPSE = 1;
224     public static final int NULL_POLICY_EXPAND = 2;
225     private int size;
226 
227     public String getField(){
228       return this.collapseField;
229     }
230 
231     public void setCache(boolean cache) {
232 
233     }
234 
235     public void setCacheSep(boolean cacheSep) {
236 
237     }
238 
239     public boolean getCacheSep() {
240       return false;
241     }
242 
243     public boolean getCache() {
244       return false;
245     }
246 
247     public int hashCode() {
248       int hashCode = super.hashCode();
249       hashCode = 31 * hashCode + collapseField.hashCode();
250       hashCode = 31 * hashCode + groupHeadSelector.hashCode();
251       hashCode = 31 * hashCode + nullPolicy;
252       return hashCode;
253     }
254 
255     public boolean equals(Object o) {
256       if (super.equals(o) == false) {
257         return false;
258       }
259 
260       if(o instanceof CollapsingPostFilter) {
261         CollapsingPostFilter c = (CollapsingPostFilter)o;
262         if(this.collapseField.equals(c.collapseField) &&
263            this.groupHeadSelector.equals(c.groupHeadSelector) &&
264            this.nullPolicy == c.nullPolicy) {
265           return true;
266         }
267       }
268       return false;
269     }
270 
271     public int getCost() {
272       return Math.max(super.getCost(), 100);
273     }
274 
275     public String toString(String s) {
276       return s;
277     }
278 
279     public CollapsingPostFilter(SolrParams localParams, SolrParams params, SolrQueryRequest request) throws IOException {
280       this.collapseField = localParams.get("field");
281       if (this.collapseField == null) {
282         throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Required 'field' param is missing.");
283       }
284 
285       this.groupHeadSelector = GroupHeadSelector.build(localParams);
286       
287       if (groupHeadSelector.type.equals(GroupHeadSelectorType.SORT) &&
288           CollapseScore.wantsCScore(groupHeadSelector.selectorText)) {
289         // we can't support Sorts that wrap functions that include "cscore()" because
290         // the abstraction layer for Sort/SortField rewriting gives each clause it's own
291         // context Map which we don't have access to -- so for now, give a useful error
292         // (as early as possible) if attempted
293         throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
294                                 "Using cscore() as a function in the 'sort' local "+
295                                 "param of the collapse parser is not supported");
296       }
297 
298         
299       
300       this.sortSpec = GroupHeadSelectorType.SORT.equals(groupHeadSelector.type) 
301         ? SortSpecParsing.parseSortSpec(groupHeadSelector.selectorText, request)
302         : null;
303 
304       this.hint = localParams.get("hint");
305       this.size = localParams.getInt("size", 100000); //Only used for collapsing on int fields.
306 
307       {
308         final SolrRequestInfo info = SolrRequestInfo.getRequestInfo();
309         assert null != info;
310         
311         // may be null in some esoteric corner usages
312         final ResponseBuilder rb = info.getResponseBuilder();
313         final SortSpec topSort = null == rb ? null : rb.getSortSpec();
314         
315         this.needsScores =
316           (info.getRsp().getReturnFields().wantsScore() ||
317            (null != topSort && topSort.includesScore()) ||
318            GroupHeadSelectorType.SCORE.equals(groupHeadSelector.type) ||
319            (GroupHeadSelectorType.SORT.equals(groupHeadSelector.type)
320             && this.sortSpec.includesScore()) ||
321            (GroupHeadSelectorType.MIN_MAX.contains(groupHeadSelector.type)
322             && CollapseScore.wantsCScore(groupHeadSelector.selectorText)) ||
323            (this.boosted != null));
324 
325         if (this.needsScores && null != rb) {
326           // regardless of why we need scores ensure the IndexSearcher will compute them
327           // for the "real" docs.  (ie: maybe we need them because we were
328           // asked to compute them for the collapsed docs, maybe we need them because in
329           // order to find the groupHead we need them computed for us.
330           
331           rb.setFieldFlags( rb.getFieldFlags() | SolrIndexSearcher.GET_SCORES);
332         }
333       }
334       
335       String nPolicy = localParams.get("nullPolicy", NULL_IGNORE);
336       if(nPolicy.equals(NULL_IGNORE)) {
337         this.nullPolicy = NULL_POLICY_IGNORE;
338       } else if (nPolicy.equals(NULL_COLLAPSE)) {
339         this.nullPolicy = NULL_POLICY_COLLAPSE;
340       } else if(nPolicy.equals((NULL_EXPAND))) {
341         this.nullPolicy = NULL_POLICY_EXPAND;
342       } else {
343         throw new IOException("Invalid nullPolicy:"+nPolicy);
344       }
345     }
346 
347     private IntIntHashMap getBoostDocs(SolrIndexSearcher indexSearcher, Map<BytesRef, Integer> boosted, Map context) throws IOException {
348       IntIntHashMap boostDocs = QueryElevationComponent.getBoostDocs(indexSearcher, boosted, context);
349       return boostDocs;
350     }
351 
352     public DelegatingCollector getFilterCollector(IndexSearcher indexSearcher) {
353       try {
354 
355         SolrIndexSearcher searcher = (SolrIndexSearcher)indexSearcher;
356         CollectorFactory collectorFactory = new CollectorFactory();
357         //Deal with boosted docs.
358         //We have to deal with it here rather then the constructor because
359         //because the QueryElevationComponent runs after the Queries are constructed.
360 
361         IntIntHashMap boostDocsMap = null;
362         Map context = null;
363         SolrRequestInfo info = SolrRequestInfo.getRequestInfo();
364         if(info != null) {
365           context = info.getReq().getContext();
366         }
367 
368         if(this.boosted == null && context != null) {
369           this.boosted = (Map<BytesRef, Integer>)context.get(QueryElevationComponent.BOOSTED_PRIORITY);
370         }
371 
372         boostDocsMap = getBoostDocs(searcher, this.boosted, context);
373         return collectorFactory.getCollector(this.collapseField,
374                                              this.groupHeadSelector,
375                                              this.sortSpec,
376                                              this.nullPolicy,
377                                              this.hint,
378                                              this.needsScores,
379                                              this.size,
380                                              boostDocsMap,
381                                              searcher);
382 
383       } catch (Exception e) {
384         throw new RuntimeException(e);
385       }
386     }
387 
388   }
389 
390   private static class ReaderWrapper extends FilterLeafReader {
391 
392     private String field;
393 
394     public ReaderWrapper(LeafReader leafReader, String field) {
395       super(leafReader);
396       this.field = field;
397     }
398 
399     public SortedDocValues getSortedDocValues(String field) {
400       return null;
401     }
402 
403     public Object getCoreCacheKey() {
404       return in.getCoreCacheKey();
405     }
406 
407     public FieldInfos getFieldInfos() {
408       Iterator<FieldInfo> it = in.getFieldInfos().iterator();
409       List<FieldInfo> newInfos = new ArrayList();
410       while(it.hasNext()) {
411         FieldInfo fieldInfo = it.next();
412 
413         if(fieldInfo.name.equals(field)) {
414           FieldInfo f = new FieldInfo(fieldInfo.name,
415                                       fieldInfo.number,
416                                       fieldInfo.hasVectors(),
417                                       fieldInfo.hasNorms(),
418                                       fieldInfo.hasPayloads(),
419                                       fieldInfo.getIndexOptions(),
420                                       DocValuesType.NONE,
421                                       fieldInfo.getDocValuesGen(),
422                                       fieldInfo.attributes());
423           newInfos.add(f);
424 
425         } else {
426           newInfos.add(fieldInfo);
427         }
428       }
429       FieldInfos infos = new FieldInfos(newInfos.toArray(new FieldInfo[newInfos.size()]));
430       return infos;
431     }
432   }
433 
434 
435   private static class DummyScorer extends Scorer {
436 
437     public float score;
438     public int docId;
439 
440     public DummyScorer() {
441       super(null);
442     }
443 
444     public float score() {
445       return score;
446     }
447 
448     public int freq() {
449       return 0;
450     }
451 
452     public int advance(int i) {
453       return -1;
454     }
455 
456     public int nextDoc() {
457       return 0;
458     }
459 
460     public int docID() {
461       return docId;
462     }
463 
464     public long cost() {
465       return 0;
466     }
467   }
468 
469 
470 
471   /*
472   * Collapses on Ordinal Values using Score to select the group head.
473   */
474 
475   private static class OrdScoreCollector extends DelegatingCollector {
476 
477     private LeafReaderContext[] contexts;
478     private FixedBitSet collapsedSet;
479     private SortedDocValues collapseValues;
480     private MultiDocValues.OrdinalMap ordinalMap;
481     private SortedDocValues segmentValues;
482     private LongValues segmentOrdinalMap;
483     private MultiDocValues.MultiSortedDocValues multiSortedDocValues;
484     private int[] ords;
485     private float[] scores;
486     private int maxDoc;
487     private int nullPolicy;
488     private float nullScore = -Float.MAX_VALUE;
489     private int nullDoc;
490     private FloatArrayList nullScores;
491     private IntArrayList boostOrds;
492     private IntArrayList boostDocs;
493     private MergeBoost mergeBoost;
494     private boolean boosts;
495 
496     public OrdScoreCollector(int maxDoc,
497                              int segments,
498                              SortedDocValues collapseValues,
499                              int nullPolicy,
500                              IntIntHashMap boostDocsMap) {
501       this.maxDoc = maxDoc;
502       this.contexts = new LeafReaderContext[segments];
503       this.collapsedSet = new FixedBitSet(maxDoc);
504       this.collapseValues = collapseValues;
505       int valueCount = collapseValues.getValueCount();
506       if(collapseValues instanceof MultiDocValues.MultiSortedDocValues) {
507         this.multiSortedDocValues = (MultiDocValues.MultiSortedDocValues)collapseValues;
508         this.ordinalMap = multiSortedDocValues.mapping;
509       }
510       this.ords = new int[valueCount];
511       Arrays.fill(this.ords, -1);
512       this.scores = new float[valueCount];
513       Arrays.fill(this.scores, -Float.MAX_VALUE);
514       this.nullPolicy = nullPolicy;
515       if(nullPolicy == CollapsingPostFilter.NULL_POLICY_EXPAND) {
516         nullScores = new FloatArrayList();
517       }
518 
519       if(boostDocsMap != null) {
520         this.boosts = true;
521         this.boostOrds = new IntArrayList();
522         this.boostDocs = new IntArrayList();
523         int[] bd = new int[boostDocsMap.size()];
524         Iterator<IntIntCursor> it =  boostDocsMap.iterator();
525         int index = -1;
526         while(it.hasNext()) {
527           IntIntCursor cursor = it.next();
528           bd[++index] = cursor.key;
529         }
530 
531         Arrays.sort(bd);
532         this.mergeBoost = new MergeBoost(bd);
533       }
534     }
535 
536     @Override
537     protected void doSetNextReader(LeafReaderContext context) throws IOException {
538       this.contexts[context.ord] = context;
539       this.docBase = context.docBase;
540       if(ordinalMap != null) {
541         this.segmentValues = this.multiSortedDocValues.values[context.ord];
542         this.segmentOrdinalMap = ordinalMap.getGlobalOrds(context.ord);
543       } else {
544         this.segmentValues = collapseValues;
545       }
546     }
547 
548     @Override
549     public void collect(int contextDoc) throws IOException {
550       int globalDoc = contextDoc+this.docBase;
551       int ord = -1;
552       if(this.ordinalMap != null) {
553         //Handle ordinalMapping case
554         ord = segmentValues.getOrd(contextDoc);
555         if(ord > -1) {
556           ord = (int)segmentOrdinalMap.get(ord);
557         }
558       } else {
559         //Handle top Level FieldCache or Single Segment Case
560         ord = segmentValues.getOrd(globalDoc);
561       }
562 
563       // Check to see if we have documents boosted by the QueryElevationComponent
564       if(boosts && mergeBoost.boost(globalDoc)) {
565         boostDocs.add(globalDoc);
566         boostOrds.add(ord);
567         return;
568       }
569 
570       if(ord > -1) {
571         float score = scorer.score();
572         if(score > scores[ord]) {
573           ords[ord] = globalDoc;
574           scores[ord] = score;
575         }
576       } else if(nullPolicy == CollapsingPostFilter.NULL_POLICY_COLLAPSE) {
577         float score = scorer.score();
578         if(score > nullScore) {
579           nullScore = score;
580           nullDoc = globalDoc;
581         }
582       } else if(nullPolicy == CollapsingPostFilter.NULL_POLICY_EXPAND) {
583         collapsedSet.set(globalDoc);
584         nullScores.add(scorer.score());
585       }
586     }
587 
588     @Override
589     public void finish() throws IOException {
590       if(contexts.length == 0) {
591         return;
592       }
593 
594       if(nullScore > 0) {
595         collapsedSet.set(nullDoc);
596       }
597 
598       //Handle the boosted docs.
599       if(this.boostOrds != null) {
600         int s = boostOrds.size();
601         for(int i=0; i<s; i++) {
602           int ord = this.boostOrds.get(i);
603           if(ord > -1) {
604             //Remove any group heads that are in the same groups as boosted documents.
605             ords[ord] = -1;
606           }
607           //Add the boosted docs to the collapsedSet
608           this.collapsedSet.set(boostDocs.get(i));
609         }
610         mergeBoost.reset(); // Reset mergeBoost because we're going to use it again.
611       }
612 
613       //Build the sorted DocSet of group heads.
614       for(int i=0; i<ords.length; i++) {
615         int doc = ords[i];
616         if(doc > -1) {
617           collapsedSet.set(doc);
618         }
619       }
620 
621       int currentContext = 0;
622       int currentDocBase = 0;
623 
624       if(ordinalMap != null) {
625         this.segmentValues = this.multiSortedDocValues.values[currentContext];
626         this.segmentOrdinalMap = this.ordinalMap.getGlobalOrds(currentContext);
627       } else {
628         this.segmentValues = collapseValues;
629       }
630 
631       int nextDocBase = currentContext+1 < contexts.length ? contexts[currentContext+1].docBase : maxDoc;
632       leafDelegate = delegate.getLeafCollector(contexts[currentContext]);
633       DummyScorer dummy = new DummyScorer();
634       leafDelegate.setScorer(dummy);
635       DocIdSetIterator it = new BitSetIterator(collapsedSet, 0L); // cost is not useful here
636       int docId = -1;
637       int index = -1;
638       while((docId = it.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
639 
640         while(docId >= nextDocBase) {
641           currentContext++;
642           currentDocBase = contexts[currentContext].docBase;
643           nextDocBase = currentContext+1 < contexts.length ? contexts[currentContext+1].docBase : maxDoc;
644           leafDelegate = delegate.getLeafCollector(contexts[currentContext]);
645           leafDelegate.setScorer(dummy);
646           if(ordinalMap != null) {
647             this.segmentValues = this.multiSortedDocValues.values[currentContext];
648             this.segmentOrdinalMap = this.ordinalMap.getGlobalOrds(currentContext);
649           }
650         }
651 
652         int contextDoc = docId-currentDocBase;
653 
654         int ord = -1;
655         if(this.ordinalMap != null) {
656           //Handle ordinalMapping case
657           ord = segmentValues.getOrd(contextDoc);
658           if(ord > -1) {
659             ord = (int)segmentOrdinalMap.get(ord);
660           }
661         } else {
662           //Handle top Level FieldCache or Single Segment Case
663           ord = segmentValues.getOrd(docId);
664         }
665 
666         if(ord > -1) {
667           dummy.score = scores[ord];
668         } else if(boosts && mergeBoost.boost(docId)) {
669           //Ignore so it doesn't mess up the null scoring.
670         } else if(this.nullPolicy == CollapsingPostFilter.NULL_POLICY_COLLAPSE) {
671           dummy.score = nullScore;
672         } else if(this.nullPolicy == CollapsingPostFilter.NULL_POLICY_EXPAND) {
673           dummy.score = nullScores.get(++index);
674         }
675 
676         dummy.docId = contextDoc;
677         leafDelegate.collect(contextDoc);
678       }
679 
680       if(delegate instanceof DelegatingCollector) {
681         ((DelegatingCollector) delegate).finish();
682       }
683     }
684   }
685 
686   /*
687   * Collapses on an integer field using the score to select the group head.
688   */
689 
690   private static class IntScoreCollector extends DelegatingCollector {
691 
692     private LeafReaderContext[] contexts;
693     private FixedBitSet collapsedSet;
694     private NumericDocValues collapseValues;
695     private IntLongHashMap cmap;
696     private int maxDoc;
697     private int nullPolicy;
698     private float nullScore = -Float.MAX_VALUE;
699     private int nullDoc;
700     private FloatArrayList nullScores;
701     private IntArrayList boostKeys;
702     private IntArrayList boostDocs;
703     private MergeBoost mergeBoost;
704     private boolean boosts;
705     private String field;
706     private int nullValue;
707 
708     public IntScoreCollector(int maxDoc,
709                              int segments,
710                              int nullValue,
711                              int nullPolicy,
712                              int size,
713                              String field,
714                              IntIntHashMap boostDocsMap) {
715       this.maxDoc = maxDoc;
716       this.contexts = new LeafReaderContext[segments];
717       this.collapsedSet = new FixedBitSet(maxDoc);
718       this.nullValue = nullValue;
719       this.nullPolicy = nullPolicy;
720       if(nullPolicy == CollapsingPostFilter.NULL_POLICY_EXPAND) {
721         nullScores = new FloatArrayList();
722       }
723       this.cmap = new IntLongHashMap(size);
724       this.field = field;
725 
726       if(boostDocsMap != null) {
727         this.boosts = true;
728         this.boostDocs = new IntArrayList();
729         this.boostKeys = new IntArrayList();
730         int[] bd = new int[boostDocsMap.size()];
731         Iterator<IntIntCursor> it =  boostDocsMap.iterator();
732         int index = -1;
733         while(it.hasNext()) {
734           IntIntCursor cursor = it.next();
735           bd[++index] = cursor.key;
736         }
737 
738         Arrays.sort(bd);
739         this.mergeBoost = new MergeBoost(bd);
740         this.boosts = true;
741       }
742 
743     }
744 
745     @Override
746     protected void doSetNextReader(LeafReaderContext context) throws IOException {
747       this.contexts[context.ord] = context;
748       this.docBase = context.docBase;
749       this.collapseValues = DocValues.getNumeric(context.reader(), this.field);
750     }
751 
752     @Override
753     public void collect(int contextDoc) throws IOException {
754 
755       int collapseValue = (int)this.collapseValues.get(contextDoc);
756       int globalDoc = docBase+contextDoc;
757 
758       // Check to see of we have documents boosted by the QueryElevationComponent
759       if(boosts && mergeBoost.boost(globalDoc)) {
760         boostDocs.add(globalDoc);
761         boostKeys.add(collapseValue);
762         return;
763       }
764 
765       if(collapseValue != nullValue) {
766         float score = scorer.score();
767         final int idx;
768         if((idx = cmap.indexOf(collapseValue)) >= 0) {
769           long scoreDoc = cmap.indexGet(idx);
770           int testScore = (int)(scoreDoc>>32);
771           int currentScore = Float.floatToRawIntBits(score);
772           if(currentScore > testScore) {
773             //Current score is higher so replace the old scoreDoc with the current scoreDoc
774             cmap.indexReplace(idx, (((long)currentScore)<<32)+globalDoc);
775           }
776         } else {
777           //Combine the score and document into a long.
778           long scoreDoc = (((long)Float.floatToRawIntBits(score))<<32)+globalDoc;
779           cmap.indexInsert(idx, collapseValue, scoreDoc);
780         }
781       } else if(nullPolicy == CollapsingPostFilter.NULL_POLICY_COLLAPSE) {
782         float score = scorer.score();
783         if(score > this.nullScore) {
784           this.nullScore = score;
785           this.nullDoc = globalDoc;
786         }
787       } else if(nullPolicy == CollapsingPostFilter.NULL_POLICY_EXPAND) {
788         collapsedSet.set(globalDoc);
789         nullScores.add(scorer.score());
790       }
791     }
792 
793     @Override
794     public void finish() throws IOException {
795       if(contexts.length == 0) {
796         return;
797       }
798 
799       if(nullScore > -1) {
800         collapsedSet.set(nullDoc);
801       }
802 
803       //Handle the boosted docs.
804       if(this.boostKeys != null) {
805         int s = boostKeys.size();
806         for(int i=0; i<s; i++) {
807           int key = this.boostKeys.get(i);
808           if(key != nullValue) {
809             cmap.remove(key);
810           }
811           //Add the boosted docs to the collapsedSet
812           this.collapsedSet.set(boostDocs.get(i));
813         }
814       }
815 
816       Iterator<IntLongCursor> it1 = cmap.iterator();
817 
818       while(it1.hasNext()) {
819         IntLongCursor cursor = it1.next();
820         int doc = (int)cursor.value;
821         collapsedSet.set(doc);
822       }
823 
824       int currentContext = 0;
825       int currentDocBase = 0;
826 
827       collapseValues = DocValues.getNumeric(contexts[currentContext].reader(), this.field);
828       int nextDocBase = currentContext+1 < contexts.length ? contexts[currentContext+1].docBase : maxDoc;
829       leafDelegate = delegate.getLeafCollector(contexts[currentContext]);
830       DummyScorer dummy = new DummyScorer();
831       leafDelegate.setScorer(dummy);
832       DocIdSetIterator it = new BitSetIterator(collapsedSet, 0L); // cost is not useful here
833       int globalDoc = -1;
834       int nullScoreIndex = 0;
835       while((globalDoc = it.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
836 
837         while(globalDoc >= nextDocBase) {
838           currentContext++;
839           currentDocBase = contexts[currentContext].docBase;
840           nextDocBase = currentContext+1 < contexts.length ? contexts[currentContext+1].docBase : maxDoc;
841           leafDelegate = delegate.getLeafCollector(contexts[currentContext]);
842           leafDelegate.setScorer(dummy);
843           collapseValues = DocValues.getNumeric(contexts[currentContext].reader(), this.field);
844         }
845 
846         int contextDoc = globalDoc-currentDocBase;
847 
848         int collapseValue = (int)collapseValues.get(contextDoc);
849         if(collapseValue != nullValue) {
850           long scoreDoc = cmap.get(collapseValue);
851           dummy.score = Float.intBitsToFloat((int)(scoreDoc>>32));
852         } else if(boosts && mergeBoost.boost(globalDoc)) {
853           //Ignore so boosted documents don't mess up the null scoring policies.
854         } else if (nullPolicy == CollapsingPostFilter.NULL_POLICY_COLLAPSE) {
855           dummy.score = nullScore;
856         } else if(nullPolicy == CollapsingPostFilter.NULL_POLICY_EXPAND) {
857           dummy.score = nullScores.get(nullScoreIndex++);
858         }
859 
860         dummy.docId = contextDoc;
861         leafDelegate.collect(contextDoc);
862       }
863 
864       if(delegate instanceof DelegatingCollector) {
865         ((DelegatingCollector) delegate).finish();
866       }
867     }
868   }
869   
870   /**
871    * Collapse on Ordinal value field.
872    */
873   private static class OrdFieldValueCollector extends DelegatingCollector {
874     private LeafReaderContext[] contexts;
875     private SortedDocValues collapseValues;
876     protected MultiDocValues.OrdinalMap ordinalMap;
877     protected SortedDocValues segmentValues;
878     protected LongValues segmentOrdinalMap;
879     protected MultiDocValues.MultiSortedDocValues multiSortedDocValues;
880 
881     private int maxDoc;
882     private int nullPolicy;
883 
884     private OrdFieldValueStrategy collapseStrategy;
885     private boolean needsScores;
886 
887     public OrdFieldValueCollector(int maxDoc,
888                                   int segments,
889                                   SortedDocValues collapseValues,
890                                   int nullPolicy,
891                                   GroupHeadSelector groupHeadSelector,
892                                   SortSpec sortSpec,
893                                   boolean needsScores,
894                                   FieldType fieldType,
895                                   IntIntHashMap boostDocs,
896                                   FunctionQuery funcQuery, IndexSearcher searcher) throws IOException{
897 
898       assert ! GroupHeadSelectorType.SCORE.equals(groupHeadSelector.type);
899       
900       this.maxDoc = maxDoc;
901       this.contexts = new LeafReaderContext[segments];
902       this.collapseValues = collapseValues;
903       if(collapseValues instanceof MultiDocValues.MultiSortedDocValues) {
904         this.multiSortedDocValues = (MultiDocValues.MultiSortedDocValues)collapseValues;
905         this.ordinalMap = multiSortedDocValues.mapping;
906       }
907 
908       int valueCount = collapseValues.getValueCount();
909       this.nullPolicy = nullPolicy;
910       this.needsScores = needsScores;
911       if (null != sortSpec) {
912         this.collapseStrategy = new OrdSortSpecStrategy(maxDoc, nullPolicy, new int[valueCount], groupHeadSelector, this.needsScores, boostDocs, sortSpec, searcher, collapseValues);
913       } else if (funcQuery != null) {
914         this.collapseStrategy =  new OrdValueSourceStrategy(maxDoc, nullPolicy, new int[valueCount], groupHeadSelector, this.needsScores, boostDocs, funcQuery, searcher, collapseValues);
915       } else {
916         if(fieldType instanceof TrieIntField) {
917           this.collapseStrategy = new OrdIntStrategy(maxDoc, nullPolicy, new int[valueCount], groupHeadSelector, this.needsScores, boostDocs, collapseValues);
918         } else if(fieldType instanceof TrieFloatField) {
919           this.collapseStrategy = new OrdFloatStrategy(maxDoc, nullPolicy, new int[valueCount], groupHeadSelector, this.needsScores, boostDocs, collapseValues);
920         } else if(fieldType instanceof TrieLongField) {
921           this.collapseStrategy =  new OrdLongStrategy(maxDoc, nullPolicy, new int[valueCount], groupHeadSelector, this.needsScores, boostDocs, collapseValues);
922         } else {
923           throw new IOException("min/max must be either TrieInt, TrieLong, TrieFloat.");
924         }
925       }
926     }
927 
928     public boolean acceptsDocsOutOfOrder() {
929       //Documents must be sent in order to this collector.
930       return false;
931     }
932 
933     public void setScorer(Scorer scorer) {
934       this.collapseStrategy.setScorer(scorer);
935     }
936 
937     public void doSetNextReader(LeafReaderContext context) throws IOException {
938       this.contexts[context.ord] = context;
939       this.docBase = context.docBase;
940       this.collapseStrategy.setNextReader(context);
941       if(ordinalMap != null) {
942         this.segmentValues = this.multiSortedDocValues.values[context.ord];
943         this.segmentOrdinalMap = ordinalMap.getGlobalOrds(context.ord);
944       } else {
945         this.segmentValues = collapseValues;
946       }
947     }
948 
949     public void collect(int contextDoc) throws IOException {
950       int globalDoc = contextDoc+this.docBase;
951       int ord = -1;
952       if(this.ordinalMap != null) {
953         ord = segmentValues.getOrd(contextDoc);
954         if(ord > -1) {
955           ord = (int)segmentOrdinalMap.get(ord);
956         }
957       } else {
958         ord = segmentValues.getOrd(globalDoc);
959       }
960       collapseStrategy.collapse(ord, contextDoc, globalDoc);
961     }
962 
963     public void finish() throws IOException {
964       if(contexts.length == 0) {
965         return;
966       }
967 
968       int currentContext = 0;
969       int currentDocBase = 0;
970 
971       if(ordinalMap != null) {
972         this.segmentValues = this.multiSortedDocValues.values[currentContext];
973         this.segmentOrdinalMap = this.ordinalMap.getGlobalOrds(currentContext);
974       } else {
975         this.segmentValues = collapseValues;
976       }
977 
978       int nextDocBase = currentContext+1 < contexts.length ? contexts[currentContext+1].docBase : maxDoc;
979       leafDelegate = delegate.getLeafCollector(contexts[currentContext]);
980       DummyScorer dummy = new DummyScorer();
981       leafDelegate.setScorer(dummy);
982       DocIdSetIterator it = new BitSetIterator(collapseStrategy.getCollapsedSet(), 0); // cost is not useful here
983       int globalDoc = -1;
984       int nullScoreIndex = 0;
985       float[] scores = collapseStrategy.getScores();
986       FloatArrayList nullScores = collapseStrategy.getNullScores();
987       float nullScore = collapseStrategy.getNullScore();
988 
989       MergeBoost mergeBoost = collapseStrategy.getMergeBoost();
990       while((globalDoc = it.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
991 
992         while(globalDoc >= nextDocBase) {
993           currentContext++;
994           currentDocBase = contexts[currentContext].docBase;
995           nextDocBase = currentContext+1 < contexts.length ? contexts[currentContext+1].docBase : maxDoc;
996           leafDelegate = delegate.getLeafCollector(contexts[currentContext]);
997           leafDelegate.setScorer(dummy);
998           if(ordinalMap != null) {
999             this.segmentValues = this.multiSortedDocValues.values[currentContext];
1000             this.segmentOrdinalMap = this.ordinalMap.getGlobalOrds(currentContext);
1001           }
1002         }
1003 
1004         int contextDoc = globalDoc-currentDocBase;
1005 
1006         if(this.needsScores){
1007           int ord = -1;
1008           if(this.ordinalMap != null) {
1009             //Handle ordinalMapping case
1010             ord = segmentValues.getOrd(contextDoc);
1011             if(ord > -1) {
1012               ord = (int)segmentOrdinalMap.get(ord);
1013             }
1014           } else {
1015             //Handle top Level FieldCache or Single Segment Case
1016             ord = segmentValues.getOrd(globalDoc);
1017           }
1018 
1019           if(ord > -1) {
1020             dummy.score = scores[ord];
1021           } else if (mergeBoost != null && mergeBoost.boost(globalDoc)) {
1022             //It's an elevated doc so no score is needed
1023             dummy.score = 0F;
1024           } else if (nullPolicy == CollapsingPostFilter.NULL_POLICY_COLLAPSE) {
1025             dummy.score = nullScore;
1026           } else if(nullPolicy == CollapsingPostFilter.NULL_POLICY_EXPAND) {
1027             dummy.score = nullScores.get(nullScoreIndex++);
1028           }
1029         }
1030 
1031         dummy.docId = contextDoc;
1032         leafDelegate.collect(contextDoc);
1033       }
1034 
1035       if(delegate instanceof DelegatingCollector) {
1036         ((DelegatingCollector) delegate).finish();
1037       }
1038     }
1039   }
1040 
1041 
1042   /**
1043    *  Collapses on an integer field.
1044    */
1045   private static class IntFieldValueCollector extends DelegatingCollector {
1046     private LeafReaderContext[] contexts;
1047     private NumericDocValues collapseValues;
1048     private int maxDoc;
1049     private int nullValue;
1050     private int nullPolicy;
1051 
1052     private IntFieldValueStrategy collapseStrategy;
1053     private boolean needsScores;
1054     private String collapseField;
1055 
1056     public IntFieldValueCollector(int maxDoc,
1057                                   int size,
1058                                   int segments,
1059                                   int nullValue,
1060                                   int nullPolicy,
1061                                   String collapseField,
1062                                   GroupHeadSelector groupHeadSelector,
1063                                   SortSpec sortSpec,
1064                                   boolean needsScores,
1065                                   FieldType fieldType,
1066                                   IntIntHashMap boostDocsMap,
1067                                   FunctionQuery funcQuery,
1068                                   IndexSearcher searcher) throws IOException{
1069 
1070       assert ! GroupHeadSelectorType.SCORE.equals(groupHeadSelector.type);
1071       
1072       this.maxDoc = maxDoc;
1073       this.contexts = new LeafReaderContext[segments];
1074       this.collapseField = collapseField;
1075       this.nullValue = nullValue;
1076       this.nullPolicy = nullPolicy;
1077       this.needsScores = needsScores;
1078       if (null != sortSpec) {
1079         this.collapseStrategy = new IntSortSpecStrategy(maxDoc, size, collapseField, nullValue, nullPolicy, groupHeadSelector, this.needsScores, boostDocsMap, sortSpec, searcher);
1080       } else if (funcQuery != null) {
1081         this.collapseStrategy =  new IntValueSourceStrategy(maxDoc, size, collapseField, nullValue, nullPolicy, groupHeadSelector, this.needsScores, boostDocsMap, funcQuery, searcher);
1082       } else {
1083         if(fieldType instanceof TrieIntField) {
1084           this.collapseStrategy = new IntIntStrategy(maxDoc, size, collapseField, nullValue, nullPolicy, groupHeadSelector, this.needsScores, boostDocsMap);
1085         } else if(fieldType instanceof TrieFloatField) {
1086           this.collapseStrategy = new IntFloatStrategy(maxDoc, size, collapseField, nullValue, nullPolicy, groupHeadSelector, this.needsScores, boostDocsMap);
1087         } else {
1088           throw new IOException("min/max must be TrieInt or TrieFloat when collapsing on numeric fields .");
1089         }
1090       }
1091     }
1092 
1093     public boolean acceptsDocsOutOfOrder() {
1094       //Documents must be sent in order to this collector.
1095       return false;
1096     }
1097 
1098     public void setScorer(Scorer scorer) {
1099       this.collapseStrategy.setScorer(scorer);
1100     }
1101 
1102     public void doSetNextReader(LeafReaderContext context) throws IOException {
1103       this.contexts[context.ord] = context;
1104       this.docBase = context.docBase;
1105       this.collapseStrategy.setNextReader(context);
1106       this.collapseValues = DocValues.getNumeric(context.reader(), this.collapseField);
1107     }
1108 
1109     public void collect(int contextDoc) throws IOException {
1110       int globalDoc = contextDoc+this.docBase;
1111       int collapseKey = (int)this.collapseValues.get(contextDoc);
1112       collapseStrategy.collapse(collapseKey, contextDoc, globalDoc);
1113     }
1114 
1115     public void finish() throws IOException {
1116       if(contexts.length == 0) {
1117         return;
1118       }
1119 
1120       int currentContext = 0;
1121       int currentDocBase = 0;
1122       this.collapseValues = DocValues.getNumeric(contexts[currentContext].reader(), this.collapseField);
1123       int nextDocBase = currentContext+1 < contexts.length ? contexts[currentContext+1].docBase : maxDoc;
1124       leafDelegate = delegate.getLeafCollector(contexts[currentContext]);
1125       DummyScorer dummy = new DummyScorer();
1126       leafDelegate.setScorer(dummy);
1127       DocIdSetIterator it = new BitSetIterator(collapseStrategy.getCollapsedSet(), 0); // cost is not useful here
1128       int globalDoc = -1;
1129       int nullScoreIndex = 0;
1130       IntIntHashMap cmap = collapseStrategy.getCollapseMap();
1131       int[] docs = collapseStrategy.getDocs();
1132       float[] scores = collapseStrategy.getScores();
1133       FloatArrayList nullScores = collapseStrategy.getNullScores();
1134       MergeBoost mergeBoost = collapseStrategy.getMergeBoost();
1135       float nullScore = collapseStrategy.getNullScore();
1136 
1137       while((globalDoc = it.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
1138 
1139         while(globalDoc >= nextDocBase) {
1140           currentContext++;
1141           currentDocBase = contexts[currentContext].docBase;
1142           nextDocBase = currentContext+1 < contexts.length ? contexts[currentContext+1].docBase : maxDoc;
1143           leafDelegate = delegate.getLeafCollector(contexts[currentContext]);
1144           leafDelegate.setScorer(dummy);
1145           this.collapseValues = DocValues.getNumeric(contexts[currentContext].reader(), this.collapseField);
1146         }
1147 
1148         int contextDoc = globalDoc-currentDocBase;
1149 
1150         if(this.needsScores){
1151           int collapseValue = (int)collapseValues.get(contextDoc);
1152           if(collapseValue != nullValue) {
1153             int pointer = cmap.get(collapseValue);
1154             dummy.score = scores[pointer];
1155           } else if (mergeBoost != null && mergeBoost.boost(globalDoc)) {
1156             //Its an elevated doc so no score is needed
1157             dummy.score = 0F;
1158           } else if (nullPolicy == CollapsingPostFilter.NULL_POLICY_COLLAPSE) {
1159             dummy.score = nullScore;
1160           } else if(nullPolicy == CollapsingPostFilter.NULL_POLICY_EXPAND) {
1161             dummy.score = nullScores.get(nullScoreIndex++);
1162           }
1163         }
1164 
1165         dummy.docId = contextDoc;
1166         leafDelegate.collect(contextDoc);
1167       }
1168 
1169       if(delegate instanceof DelegatingCollector) {
1170         ((DelegatingCollector) delegate).finish();
1171       }
1172     }
1173   }
1174 
1175   private static class CollectorFactory {
1176 
1177     public DelegatingCollector getCollector(String collapseField,
1178                                             GroupHeadSelector groupHeadSelector,
1179                                             SortSpec sortSpec,
1180                                             int nullPolicy,
1181                                             String hint,
1182                                             boolean needsScores,
1183                                             int size,
1184                                             IntIntHashMap boostDocs,
1185                                             SolrIndexSearcher searcher) throws IOException {
1186 
1187       SortedDocValues docValues = null;
1188       FunctionQuery funcQuery = null;
1189 
1190       FieldType collapseFieldType = searcher.getSchema().getField(collapseField).getType();
1191       String defaultValue = searcher.getSchema().getField(collapseField).getDefaultValue();
1192 
1193       if(collapseFieldType instanceof StrField) {
1194         if(HINT_TOP_FC.equals(hint)) {
1195 
1196             /*
1197             * This hint forces the use of the top level field cache for String fields.
1198             * This is VERY fast at query time but slower to warm and causes insanity.
1199             */
1200 
1201           Map<String, UninvertingReader.Type> mapping = new HashMap();
1202           mapping.put(collapseField, UninvertingReader.Type.SORTED);
1203           UninvertingReader uninvertingReader = new UninvertingReader(new ReaderWrapper(searcher.getLeafReader(), collapseField), mapping);
1204           docValues = uninvertingReader.getSortedDocValues(collapseField);
1205         } else {
1206           docValues = DocValues.getSorted(searcher.getLeafReader(), collapseField);
1207         }
1208       } else {
1209         if(HINT_TOP_FC.equals(hint)) {
1210           throw new IOException("top_fc hint is only supported when collapsing on String Fields");
1211         }
1212       }
1213 
1214       FieldType minMaxFieldType = null;
1215       if (GroupHeadSelectorType.MIN_MAX.contains(groupHeadSelector.type)) {
1216         final String text = groupHeadSelector.selectorText;
1217         if (text.indexOf("(") == -1) {
1218           minMaxFieldType = searcher.getSchema().getField(text).getType();
1219         } else {
1220           LocalSolrQueryRequest request = null;
1221           try {
1222             SolrParams params = new ModifiableSolrParams();
1223             request = new LocalSolrQueryRequest(searcher.getCore(), params);
1224             FunctionQParser functionQParser = new FunctionQParser(text, null, null,request);
1225             funcQuery = (FunctionQuery)functionQParser.parse();
1226           } catch (Exception e) {
1227             throw new IOException(e);
1228           } finally {
1229             request.close();
1230           }
1231         }
1232       }
1233 
1234       int maxDoc = searcher.maxDoc();
1235       int leafCount = searcher.getTopReaderContext().leaves().size();
1236 
1237       if (GroupHeadSelectorType.SCORE.equals(groupHeadSelector.type)) {
1238         
1239         if (collapseFieldType instanceof StrField) {
1240 
1241           return new OrdScoreCollector(maxDoc, leafCount, docValues, nullPolicy, boostDocs);
1242 
1243         } else if (collapseFieldType instanceof TrieIntField ||
1244                    collapseFieldType instanceof TrieFloatField) {
1245 
1246           int nullValue = 0;
1247 
1248           if(collapseFieldType instanceof TrieFloatField) {
1249             if(defaultValue != null) {
1250               nullValue = Float.floatToIntBits(Float.parseFloat(defaultValue));
1251             } else {
1252               nullValue = Float.floatToIntBits(0.0f);
1253             }
1254           } else {
1255             if(defaultValue != null) {
1256               nullValue = Integer.parseInt(defaultValue);
1257             }
1258           }
1259 
1260           return new IntScoreCollector(maxDoc, leafCount, nullValue, nullPolicy, size, collapseField, boostDocs);
1261 
1262         } else {
1263           throw new IOException("64 bit numeric collapse fields are not supported");
1264         }
1265         
1266       } else { // min, max, sort, etc.. something other then just "score"
1267 
1268         if (collapseFieldType instanceof StrField) {
1269 
1270           return new OrdFieldValueCollector(maxDoc,
1271                                             leafCount,
1272                                             docValues,
1273                                             nullPolicy,
1274                                             groupHeadSelector,
1275                                             sortSpec,
1276                                             needsScores,
1277                                             minMaxFieldType,
1278                                             boostDocs,
1279                                             funcQuery,
1280                                             searcher);
1281 
1282         } else if((collapseFieldType instanceof TrieIntField ||
1283                    collapseFieldType instanceof TrieFloatField)) {
1284 
1285           int nullValue = 0;
1286 
1287           if(collapseFieldType instanceof TrieFloatField) {
1288             if(defaultValue != null) {
1289               nullValue = Float.floatToIntBits(Float.parseFloat(defaultValue));
1290             } else {
1291               nullValue = Float.floatToIntBits(0.0f);
1292             }
1293           } else {
1294             if(defaultValue != null) {
1295               nullValue = Integer.parseInt(defaultValue);
1296             }
1297           }
1298 
1299           return new IntFieldValueCollector(maxDoc,
1300                                             size,
1301                                             leafCount,
1302                                             nullValue,
1303                                             nullPolicy,
1304                                             collapseField,
1305                                             groupHeadSelector,
1306                                             sortSpec,
1307                                             needsScores,
1308                                             minMaxFieldType,
1309                                             boostDocs,
1310                                             funcQuery,
1311                                             searcher);
1312         } else {
1313           throw new IOException("64 bit numeric collapse fields are not supported");
1314         }
1315         
1316       }
1317     }
1318   }
1319 
1320   public static final class CollapseScore {
1321     /** 
1322      * Inspects the GroupHeadSelector to determine if this CollapseScore is needed.
1323      * If it is, then "this" will be added to the readerContext 
1324      * using the "CSCORE" key, and true will be returned.  If not returns false.
1325      */
1326     public boolean setupIfNeeded(final GroupHeadSelector groupHeadSelector,
1327                                  final Map readerContext) {
1328       // HACK, but not really any better options until/unless we can recursively
1329       // ask value sources if they depend on score
1330       if (wantsCScore(groupHeadSelector.selectorText)) {
1331         readerContext.put("CSCORE", this);
1332         return true;
1333       }
1334       return false;
1335     }
1336 
1337     /** 
1338      * Huge HACK, but not really any better options until/unless we can recursively 
1339      * ask value sources if they depend on score 
1340      */
1341     public static boolean wantsCScore(final String text) {
1342       return (0 <= text.indexOf("cscore()"));
1343     }
1344     
1345     private CollapseScore() {
1346       // No-Op
1347     }
1348     
1349     public float score;
1350   }
1351 
1352 
1353   /*
1354   * Collapse Strategies
1355   */
1356 
1357   /**
1358    * The abstract base Strategy for collapse strategies that collapse on an ordinal
1359    * using min/max field value to select the group head.
1360    *
1361    */
1362   private static abstract class OrdFieldValueStrategy {
1363     protected int nullPolicy;
1364     protected int[] ords; 
1365     protected Scorer scorer;
1366     protected FloatArrayList nullScores;
1367     protected float nullScore;
1368     protected float[] scores;
1369     protected FixedBitSet collapsedSet;
1370     protected int nullDoc = -1;
1371     protected boolean needsScores;
1372     protected boolean boosts;
1373     protected IntArrayList boostOrds;
1374     protected IntArrayList boostDocs;
1375     protected MergeBoost mergeBoost;
1376     protected boolean boosted;
1377 
1378     public abstract void collapse(int ord, int contextDoc, int globalDoc) throws IOException;
1379     public abstract void setNextReader(LeafReaderContext context) throws IOException;
1380 
1381     public OrdFieldValueStrategy(int maxDoc,
1382                                  int[] ords,
1383                                  int nullPolicy,
1384                                  boolean needsScores,
1385                                  IntIntHashMap boostDocsMap,
1386                                  SortedDocValues values) {
1387       this.ords = ords;
1388       Arrays.fill(ords, -1);
1389       this.nullPolicy = nullPolicy;
1390       this.needsScores = needsScores;
1391       this.collapsedSet = new FixedBitSet(maxDoc);
1392       if(boostDocsMap != null) {
1393         this.boosts = true;
1394         this.boostOrds = new IntArrayList();
1395         this.boostDocs = new IntArrayList();
1396         int[] bd = new int[boostDocsMap.size()];
1397         Iterator<IntIntCursor> it =  boostDocsMap.iterator();
1398         int index = -1;
1399         while(it.hasNext()) {
1400           IntIntCursor cursor = it.next();
1401           bd[++index] = cursor.key;
1402         }
1403 
1404         Arrays.sort(bd);
1405         this.mergeBoost = new MergeBoost(bd);
1406         this.boosted = true;
1407       }
1408     }
1409 
1410     public MergeBoost getMergeBoost() {
1411       return this.mergeBoost;
1412     }
1413 
1414     public FixedBitSet getCollapsedSet() {
1415       if(nullDoc > -1) {
1416         this.collapsedSet.set(nullDoc);
1417       }
1418 
1419       if(this.boostOrds != null) {
1420         int s = boostOrds.size();
1421         for(int i=0; i<s; i++) {
1422           int ord = boostOrds.get(i);
1423           if(ord > -1) {
1424             ords[ord] = -1;
1425           }
1426           collapsedSet.set(boostDocs.get(i));
1427         }
1428 
1429         mergeBoost.reset();
1430       }
1431 
1432       for(int i=0; i<ords.length; i++) {
1433         int doc = ords[i];
1434         if(doc > -1) {
1435           collapsedSet.set(doc);
1436         }
1437       }
1438 
1439       return collapsedSet;
1440     }
1441 
1442     public void setScorer(Scorer scorer) {
1443       this.scorer = scorer;
1444     }
1445 
1446     public FloatArrayList getNullScores() {
1447       return nullScores;
1448     }
1449 
1450     public float getNullScore() {
1451       return this.nullScore;
1452     }
1453 
1454     public float[] getScores() {
1455       return scores;
1456     }
1457   }
1458 
1459   /*
1460    * Strategy for collapsing on ordinal using min/max of an int field to select the group head.
1461    */
1462   private static class OrdIntStrategy extends OrdFieldValueStrategy {
1463 
1464     private final String field;
1465     private NumericDocValues minMaxValues;
1466     private IntCompare comp;
1467     private int nullVal;
1468     private int[] ordVals;
1469 
1470     public OrdIntStrategy(int maxDoc,
1471                           int nullPolicy,
1472                           int[] ords,
1473                           GroupHeadSelector groupHeadSelector,
1474                           boolean needsScores,
1475                           IntIntHashMap boostDocs,
1476                           SortedDocValues values) throws IOException {
1477       super(maxDoc, ords, nullPolicy, needsScores, boostDocs, values);
1478       this.field = groupHeadSelector.selectorText;
1479       this.ordVals = new int[ords.length];
1480 
1481       assert GroupHeadSelectorType.MIN_MAX.contains(groupHeadSelector.type);
1482       
1483       if (GroupHeadSelectorType.MAX.equals(groupHeadSelector.type)) {
1484         comp = new MaxIntComp();
1485         Arrays.fill(ordVals, Integer.MIN_VALUE);
1486       } else {
1487         comp = new MinIntComp();
1488         Arrays.fill(ordVals, Integer.MAX_VALUE);
1489         this.nullVal = Integer.MAX_VALUE;
1490       }
1491 
1492       if(needsScores) {
1493         this.scores = new float[ords.length];
1494         if(nullPolicy == CollapsingPostFilter.NULL_POLICY_EXPAND) {
1495           nullScores = new FloatArrayList();
1496         }
1497       }
1498     }
1499 
1500     public void setNextReader(LeafReaderContext context) throws IOException {
1501       this.minMaxValues = DocValues.getNumeric(context.reader(), this.field);
1502     }
1503 
1504     public void collapse(int ord, int contextDoc, int globalDoc) throws IOException {
1505 
1506       if(this.boosted && mergeBoost.boost(globalDoc)) {
1507         this.boostDocs.add(globalDoc);
1508         this.boostOrds.add(ord);
1509         return;
1510       }
1511 
1512       int currentVal = (int) minMaxValues.get(contextDoc);
1513 
1514       if(ord > -1) {
1515         if(comp.test(currentVal, ordVals[ord])) {
1516           ords[ord] = globalDoc;
1517           ordVals[ord] = currentVal;
1518           if(needsScores) {
1519             scores[ord] = scorer.score();
1520           }
1521         }
1522       } else if(this.nullPolicy == CollapsingPostFilter.NULL_POLICY_COLLAPSE) {
1523         if(comp.test(currentVal, nullVal)) {
1524           nullVal = currentVal;
1525           nullDoc = globalDoc;
1526           if(needsScores) {
1527             nullScore = scorer.score();
1528           }
1529         }
1530       } else if(this.nullPolicy == CollapsingPostFilter.NULL_POLICY_EXPAND) {
1531         this.collapsedSet.set(globalDoc);
1532         if(needsScores) {
1533           nullScores.add(scorer.score());
1534         }
1535       }
1536     }
1537   }
1538 
1539   /**
1540    * Strategy for collapsing on ordinal and using the min/max value of a float
1541    * field to select the group head
1542    */
1543   private static class OrdFloatStrategy extends OrdFieldValueStrategy {
1544 
1545     private final String field;
1546     private NumericDocValues minMaxValues;
1547     private FloatCompare comp;
1548     private float nullVal;
1549     private float[] ordVals;
1550 
1551     public OrdFloatStrategy(int maxDoc,
1552                             int nullPolicy,
1553                             int[] ords,
1554                             GroupHeadSelector groupHeadSelector,
1555                             boolean needsScores,
1556                             IntIntHashMap boostDocs,
1557                             SortedDocValues values) throws IOException {
1558       super(maxDoc, ords, nullPolicy, needsScores, boostDocs, values);
1559       this.field = groupHeadSelector.selectorText;
1560       this.ordVals = new float[ords.length];
1561       
1562       assert GroupHeadSelectorType.MIN_MAX.contains(groupHeadSelector.type);
1563 
1564       if (GroupHeadSelectorType.MAX.equals(groupHeadSelector.type)) {
1565         comp = new MaxFloatComp();
1566         Arrays.fill(ordVals, -Float.MAX_VALUE);
1567         this.nullVal = -Float.MAX_VALUE;
1568       } else {
1569         comp = new MinFloatComp();
1570         Arrays.fill(ordVals, Float.MAX_VALUE);
1571         this.nullVal = Float.MAX_VALUE;
1572       }
1573 
1574       if(needsScores) {
1575         this.scores = new float[ords.length];
1576         if(nullPolicy == CollapsingPostFilter.NULL_POLICY_EXPAND) {
1577           nullScores = new FloatArrayList();
1578         }
1579       }
1580     }
1581 
1582     public void setNextReader(LeafReaderContext context) throws IOException {
1583       this.minMaxValues = DocValues.getNumeric(context.reader(), this.field);
1584     }
1585 
1586     public void collapse(int ord, int contextDoc, int globalDoc) throws IOException {
1587 
1588       if(this.boosted && mergeBoost.boost(globalDoc)) {
1589         this.boostDocs.add(globalDoc);
1590         this.boostOrds.add(ord);
1591         return;
1592       }
1593 
1594       int currentMinMax = (int) minMaxValues.get(contextDoc);
1595       float currentVal = Float.intBitsToFloat(currentMinMax);
1596 
1597       if(ord > -1) {
1598         if(comp.test(currentVal, ordVals[ord])) {
1599           ords[ord] = globalDoc;
1600           ordVals[ord] = currentVal;
1601           if(needsScores) {
1602             scores[ord] = scorer.score();
1603           }
1604         }
1605       } else if(this.nullPolicy == CollapsingPostFilter.NULL_POLICY_COLLAPSE) {
1606         if(comp.test(currentVal, nullVal)) {
1607           nullVal = currentVal;
1608           nullDoc = globalDoc;
1609           if(needsScores) {
1610             nullScore = scorer.score();
1611           }
1612         }
1613       } else if(this.nullPolicy == CollapsingPostFilter.NULL_POLICY_EXPAND) {
1614         this.collapsedSet.set(globalDoc);
1615         if(needsScores) {
1616           nullScores.add(scorer.score());
1617         }
1618       }
1619     }
1620   }
1621 
1622   /*
1623   * Strategy for collapsing on ordinal and using the min/max value of a long
1624   * field to select the group head
1625   */
1626 
1627   private static class OrdLongStrategy extends OrdFieldValueStrategy {
1628 
1629     private final String field;
1630     private NumericDocValues minMaxVals;
1631     private LongCompare comp;
1632     private long nullVal;
1633     private long[] ordVals;
1634 
1635     public OrdLongStrategy(int maxDoc,
1636                            int nullPolicy,
1637                            int[] ords,
1638                            GroupHeadSelector groupHeadSelector,
1639                            boolean needsScores,
1640                            IntIntHashMap boostDocs, SortedDocValues values) throws IOException {
1641       super(maxDoc, ords, nullPolicy, needsScores, boostDocs, values);
1642       this.field = groupHeadSelector.selectorText;
1643       this.ordVals = new long[ords.length];
1644 
1645       assert GroupHeadSelectorType.MIN_MAX.contains(groupHeadSelector.type);
1646       
1647       if (GroupHeadSelectorType.MAX.equals(groupHeadSelector.type)) {
1648         comp = new MaxLongComp();
1649         Arrays.fill(ordVals, Long.MIN_VALUE);
1650       } else {
1651         this.nullVal = Long.MAX_VALUE;
1652         comp = new MinLongComp();
1653         Arrays.fill(ordVals, Long.MAX_VALUE);
1654       }
1655 
1656       if(needsScores) {
1657         this.scores = new float[ords.length];
1658         if(nullPolicy == CollapsingPostFilter.NULL_POLICY_EXPAND) {
1659           nullScores = new FloatArrayList();
1660         }
1661       }
1662     }
1663 
1664     public void setNextReader(LeafReaderContext context) throws IOException {
1665       this.minMaxVals = DocValues.getNumeric(context.reader(), this.field);
1666     }
1667 
1668     public void collapse(int ord, int contextDoc, int globalDoc) throws IOException {
1669 
1670       if(boosted && mergeBoost.boost(globalDoc)) {
1671         this.boostOrds.add(ord);
1672         this.boostDocs.add(globalDoc);
1673         return;
1674       }
1675 
1676       long currentVal = minMaxVals.get(contextDoc);
1677       if(ord > -1) {
1678         if(comp.test(currentVal, ordVals[ord])) {
1679           ords[ord] = globalDoc;
1680           ordVals[ord] = currentVal;
1681           if(needsScores) {
1682             scores[ord] = scorer.score();
1683           }
1684         }
1685       } else if(this.nullPolicy == CollapsingPostFilter.NULL_POLICY_COLLAPSE) {
1686         if(comp.test(currentVal, nullVal)) {
1687           nullVal = currentVal;
1688           nullDoc = globalDoc;
1689           if(needsScores) {
1690             nullScore = scorer.score();
1691           }
1692         }
1693       } else if(this.nullPolicy == CollapsingPostFilter.NULL_POLICY_EXPAND) {
1694         this.collapsedSet.set(globalDoc);
1695         if(needsScores) {
1696           nullScores.add(scorer.score());
1697         }
1698       }
1699     }
1700   }
1701 
1702   /*
1703    * Strategy for collapsing on ordinal and using the min/max value of a value source function
1704    * to select the group head
1705    */
1706   private static class OrdValueSourceStrategy extends OrdFieldValueStrategy {
1707 
1708     private FloatCompare comp;
1709     private float nullVal;
1710     private ValueSource valueSource;
1711     private FunctionValues functionValues;
1712     private float[] ordVals;
1713     private Map rcontext;
1714     private final CollapseScore collapseScore = new CollapseScore();
1715     private final boolean cscore;
1716     private float score;
1717 
1718     public OrdValueSourceStrategy(int maxDoc,
1719                                   int nullPolicy,
1720                                   int[] ords,
1721                                   GroupHeadSelector groupHeadSelector,
1722                                   boolean needsScores,
1723                                   IntIntHashMap boostDocs,
1724                                   FunctionQuery funcQuery,
1725                                   IndexSearcher searcher,
1726                                   SortedDocValues values) throws IOException {
1727       super(maxDoc, ords, nullPolicy, needsScores, boostDocs, values);
1728       this.valueSource = funcQuery.getValueSource();
1729       this.rcontext = ValueSource.newContext(searcher);
1730       this.ordVals = new float[ords.length];
1731 
1732       assert GroupHeadSelectorType.MIN_MAX.contains(groupHeadSelector.type);
1733       
1734       if (GroupHeadSelectorType.MAX.equals(groupHeadSelector.type)) {
1735         comp = new MaxFloatComp();
1736         Arrays.fill(ordVals, -Float.MAX_VALUE );
1737       } else {
1738         this.nullVal = Float.MAX_VALUE;
1739         comp = new MinFloatComp();
1740         Arrays.fill(ordVals, Float.MAX_VALUE);
1741       }
1742 
1743       this.cscore = collapseScore.setupIfNeeded(groupHeadSelector, rcontext);
1744 
1745       if(this.needsScores) {
1746         this.scores = new float[ords.length];
1747         if(nullPolicy == CollapsingPostFilter.NULL_POLICY_EXPAND) {
1748           nullScores = new FloatArrayList();
1749         }
1750       }
1751     }
1752 
1753     public void setNextReader(LeafReaderContext context) throws IOException {
1754       functionValues = this.valueSource.getValues(rcontext, context);
1755     }
1756 
1757     public void collapse(int ord, int contextDoc, int globalDoc) throws IOException {
1758 
1759       if(boosted && mergeBoost.boost(globalDoc)) {
1760         this.boostOrds.add(ord);
1761         this.boostDocs.add(globalDoc);
1762       }
1763 
1764       if(needsScores || cscore) {
1765         this.score = scorer.score();
1766         this.collapseScore.score = score;
1767       }
1768 
1769       float currentVal = functionValues.floatVal(contextDoc);
1770 
1771       if(ord > -1) {
1772         if(comp.test(currentVal, ordVals[ord])) {
1773           ords[ord] = globalDoc;
1774           ordVals[ord] = currentVal;
1775           if(needsScores) {
1776             scores[ord] = score;
1777           }
1778         }
1779       } else if(this.nullPolicy == CollapsingPostFilter.NULL_POLICY_COLLAPSE) {
1780         if(comp.test(currentVal, nullVal)) {
1781           nullVal = currentVal;
1782           nullDoc = globalDoc;
1783           if(needsScores) {
1784             nullScore = score;
1785           }
1786         }
1787       } else if(this.nullPolicy == CollapsingPostFilter.NULL_POLICY_EXPAND) {
1788         this.collapsedSet.set(globalDoc);
1789         if(needsScores) {
1790           nullScores.add(score);
1791         }
1792       }
1793     }
1794   }
1795 
1796   /*
1797    * Strategy for collapsing on ordinal and using the first document according to a complex sort
1798    * as the group head
1799    */
1800   private static class OrdSortSpecStrategy extends OrdFieldValueStrategy {
1801 
1802     private final SortFieldsCompare compareState;
1803     private final SortSpec sortSpec;
1804     private final Sort sort;
1805 
1806     private float score;
1807 
1808     public OrdSortSpecStrategy(int maxDoc,
1809                                int nullPolicy,
1810                                int[] ords,
1811                                GroupHeadSelector groupHeadSelector,
1812                                boolean needsScores,
1813                                IntIntHashMap boostDocs,
1814                                SortSpec sortSpec,
1815                                IndexSearcher searcher,
1816                                SortedDocValues values) throws IOException {
1817       super(maxDoc, ords, nullPolicy, needsScores, boostDocs, values);
1818       
1819       assert GroupHeadSelectorType.SORT.equals(groupHeadSelector.type);
1820       
1821       this.sortSpec = sortSpec;
1822       this.sort = rewriteSort(sortSpec, searcher);
1823       
1824       this.compareState = new SortFieldsCompare(sort.getSort(), ords.length);
1825 
1826       if (this.needsScores) {
1827         this.scores = new float[ords.length];
1828         if(nullPolicy == CollapsingPostFilter.NULL_POLICY_EXPAND) {
1829           nullScores = new FloatArrayList();
1830         }
1831       }
1832     }
1833 
1834     @Override
1835     public void setNextReader(LeafReaderContext context) throws IOException {
1836       compareState.setNextReader(context);
1837     }
1838 
1839     @Override
1840     public void setScorer(Scorer s) {
1841       super.setScorer(s);
1842       this.compareState.setScorer(s);
1843     }
1844     
1845     @Override
1846     public void collapse(int ord, int contextDoc, int globalDoc) throws IOException {
1847       
1848       if(boosted && mergeBoost.boost(globalDoc)) {
1849         this.boostOrds.add(ord);
1850         this.boostDocs.add(globalDoc);
1851       }
1852 
1853       if (needsScores) {
1854         this.score = scorer.score();
1855       }
1856 
1857       if (ord > -1) { // real collapseKey
1858         if (-1 == ords[ord]) {
1859           // we've never seen this ord (aka: collapseKey) before, treat it as group head for now
1860           compareState.setGroupValues(ord, contextDoc);
1861           ords[ord] = globalDoc;
1862           if (needsScores) {
1863             scores[ord] = score;
1864           }
1865         } else {
1866           // test this ord to see if it's a new group leader
1867           if (compareState.testAndSetGroupValues(ord, contextDoc)) {
1868             ords[ord] = globalDoc;
1869             if (needsScores) {
1870               scores[ord] = score;
1871             }
1872           }
1873         }
1874       } else if (this.nullPolicy == CollapsingPostFilter.NULL_POLICY_COLLAPSE) {
1875         if (-1 == nullDoc) {
1876           // we've never seen a doc with null collapse key yet, treat it as the null group head for now
1877           compareState.setNullGroupValues(contextDoc);
1878           nullDoc = globalDoc;
1879           if (needsScores) {
1880             nullScore = score;
1881           }
1882         } else {
1883           // test this doc to see if it's the new null leader
1884           if (compareState.testAndSetNullGroupValues(contextDoc)) {
1885             nullDoc = globalDoc;
1886             if (needsScores) {
1887               nullScore = score;
1888             }
1889           }
1890         }
1891       } else if(this.nullPolicy == CollapsingPostFilter.NULL_POLICY_EXPAND) {
1892         this.collapsedSet.set(globalDoc);
1893         if (needsScores) {
1894           nullScores.add(score);
1895         }
1896       }
1897     }
1898   }
1899 
1900   /*
1901   * Base strategy for collapsing on a 32 bit numeric field and selecting a group head
1902   * based on min/max value of a 32 bit numeric field.
1903   */
1904 
1905   private static abstract class IntFieldValueStrategy {
1906     protected int nullPolicy;
1907     protected IntIntHashMap cmap;
1908     protected Scorer scorer;
1909     protected FloatArrayList nullScores;
1910     protected float nullScore;
1911     protected float[] scores;
1912     protected FixedBitSet collapsedSet;
1913     protected int nullDoc = -1;
1914     protected boolean needsScores;
1915     protected String collapseField;
1916     protected int[] docs;
1917     protected int nullValue;
1918     protected IntArrayList boostDocs;
1919     protected IntArrayList boostKeys;
1920     protected boolean boosts;
1921     protected MergeBoost mergeBoost;
1922 
1923     public abstract void collapse(int collapseKey, int contextDoc, int globalDoc) throws IOException;
1924     public abstract void setNextReader(LeafReaderContext context) throws IOException;
1925 
1926     public IntFieldValueStrategy(int maxDoc,
1927                                  int size,
1928                                  String collapseField,
1929                                  int nullValue,
1930                                  int nullPolicy,
1931                                  boolean needsScores,
1932                                  IntIntHashMap boostDocsMap) {
1933       this.collapseField = collapseField;
1934       this.nullValue = nullValue;
1935       this.nullPolicy = nullPolicy;
1936       this.needsScores = needsScores;
1937       this.collapsedSet = new FixedBitSet(maxDoc);
1938       this.cmap = new IntIntHashMap(size);
1939       this.docs = new int[size];
1940       if(boostDocsMap != null) {
1941         this.boosts = true;
1942         this.boostDocs = new IntArrayList();
1943         this.boostKeys = new IntArrayList();
1944         int[] bd = new int[boostDocsMap.size()];
1945         Iterator<IntIntCursor> it =  boostDocsMap.iterator();
1946         int index = -1;
1947         while(it.hasNext()) {
1948           IntIntCursor cursor = it.next();
1949           bd[++index] = cursor.key;
1950         }
1951 
1952         Arrays.sort(bd);
1953         this.mergeBoost = new MergeBoost(bd);
1954       }
1955     }
1956 
1957     public FixedBitSet getCollapsedSet() {
1958 
1959       if(nullDoc > -1) {
1960         this.collapsedSet.set(nullDoc);
1961       }
1962 
1963       //Handle the boosted docs.
1964       if(this.boostKeys != null) {
1965         int s = boostKeys.size();
1966         for(int i=0; i<s; i++) {
1967           int key = this.boostKeys.get(i);
1968           if(key != nullValue) {
1969             cmap.remove(key);
1970           }
1971           //Add the boosted docs to the collapsedSet
1972           this.collapsedSet.set(boostDocs.get(i));
1973         }
1974 
1975         mergeBoost.reset();
1976       }
1977 
1978       Iterator<IntIntCursor> it1 = cmap.iterator();
1979       while(it1.hasNext()) {
1980         IntIntCursor cursor = it1.next();
1981         int pointer = cursor.value;
1982         collapsedSet.set(docs[pointer]);
1983       }
1984 
1985       return collapsedSet;
1986     }
1987 
1988     public void setScorer(Scorer scorer) {
1989       this.scorer = scorer;
1990     }
1991 
1992     public FloatArrayList getNullScores() {
1993       return nullScores;
1994     }
1995 
1996     public IntIntHashMap getCollapseMap() {
1997       return cmap;
1998     }
1999 
2000     public float getNullScore() {
2001       return this.nullScore;
2002     }
2003 
2004     public float[] getScores() {
2005       return scores;
2006     }
2007 
2008     public int[] getDocs() { return docs;}
2009 
2010     public MergeBoost getMergeBoost()  {
2011       return this.mergeBoost;
2012     }
2013   }
2014 
2015   /*
2016    *  Strategy for collapsing on a 32 bit numeric field and selecting the group head based
2017    *  on the min/max value of a 32 bit field numeric field.
2018    */
2019   private static class IntIntStrategy extends IntFieldValueStrategy {
2020 
2021     private final String field;
2022     private NumericDocValues minMaxVals;
2023     private int[] testValues;
2024     private IntCompare comp;
2025     private int nullCompVal;
2026 
2027     private int index=-1;
2028 
2029     public IntIntStrategy(int maxDoc,
2030                           int size,
2031                           String collapseField,
2032                           int nullValue,
2033                           int nullPolicy,
2034                           GroupHeadSelector groupHeadSelector,
2035                           boolean needsScores,
2036                           IntIntHashMap boostDocs) throws IOException {
2037 
2038       super(maxDoc, size, collapseField, nullValue, nullPolicy, needsScores, boostDocs);
2039       this.field = groupHeadSelector.selectorText;
2040       this.testValues = new int[size];
2041 
2042       assert GroupHeadSelectorType.MIN_MAX.contains(groupHeadSelector.type);
2043       
2044       if (GroupHeadSelectorType.MAX.equals(groupHeadSelector.type)) {
2045         comp = new MaxIntComp();
2046         this.nullCompVal = Integer.MIN_VALUE;
2047       } else {
2048         comp = new MinIntComp();
2049         this.nullCompVal = Integer.MAX_VALUE;
2050       }
2051 
2052       if(needsScores) {
2053         this.scores = new float[size];
2054         if(nullPolicy == CollapsingPostFilter.NULL_POLICY_EXPAND) {
2055           nullScores = new FloatArrayList();
2056         }
2057       }
2058     }
2059 
2060     public void setNextReader(LeafReaderContext context) throws IOException {
2061       this.minMaxVals = DocValues.getNumeric(context.reader(), this.field);
2062     }
2063 
2064     public void collapse(int collapseKey, int contextDoc, int globalDoc) throws IOException {
2065 
2066       // Check to see if we have documents boosted by the QueryElevationComponent
2067       if(boosts && mergeBoost.boost(globalDoc)) {
2068         boostDocs.add(globalDoc);
2069         boostKeys.add(collapseKey);
2070         return;
2071       }
2072 
2073       int currentVal = (int) minMaxVals.get(contextDoc);
2074 
2075       if(collapseKey != nullValue) {
2076         final int idx;
2077         if((idx = cmap.indexOf(collapseKey)) >= 0) {
2078           int pointer = cmap.indexGet(idx);
2079           if(comp.test(currentVal, testValues[pointer])) {
2080             testValues[pointer]= currentVal;
2081             docs[pointer] = globalDoc;
2082             if(needsScores) {
2083               scores[pointer] = scorer.score();
2084             }
2085           }
2086         } else {
2087           ++index;
2088           cmap.put(collapseKey, index);
2089           if(index == testValues.length) {
2090             testValues = ArrayUtil.grow(testValues);
2091             docs = ArrayUtil.grow(docs);
2092             if(needsScores) {
2093               scores = ArrayUtil.grow(scores);
2094             }
2095           }
2096 
2097           testValues[index] = currentVal;
2098           docs[index] = (globalDoc);
2099 
2100           if(needsScores) {
2101             scores[index] = scorer.score();
2102           }
2103         }
2104       } else if(this.nullPolicy == CollapsingPostFilter.NULL_POLICY_COLLAPSE) {
2105         if(comp.test(currentVal, nullCompVal)) {
2106           nullCompVal = currentVal;
2107           nullDoc = globalDoc;
2108           if(needsScores) {
2109             nullScore = scorer.score();
2110           }
2111         }
2112       } else if(this.nullPolicy == CollapsingPostFilter.NULL_POLICY_EXPAND) {
2113         this.collapsedSet.set(globalDoc);
2114         if(needsScores) {
2115           nullScores.add(scorer.score());
2116         }
2117       }
2118     }
2119   }
2120 
2121   private static class IntFloatStrategy extends IntFieldValueStrategy {
2122 
2123     private final String field;
2124     private NumericDocValues minMaxVals;
2125     private float[] testValues;
2126     private FloatCompare comp;
2127     private float nullCompVal;
2128 
2129     private int index=-1;
2130 
2131     public IntFloatStrategy(int maxDoc,
2132                             int size,
2133                             String collapseField,
2134                             int nullValue,
2135                             int nullPolicy,
2136                             GroupHeadSelector groupHeadSelector,
2137                             boolean needsScores,
2138                             IntIntHashMap boostDocs) throws IOException {
2139 
2140       super(maxDoc, size, collapseField, nullValue, nullPolicy, needsScores, boostDocs);
2141       this.field = groupHeadSelector.selectorText;
2142       this.testValues = new float[size];
2143 
2144       assert GroupHeadSelectorType.MIN_MAX.contains(groupHeadSelector.type);
2145       
2146       if (GroupHeadSelectorType.MAX.equals(groupHeadSelector.type)) {
2147         comp = new MaxFloatComp();
2148         this.nullCompVal = -Float.MAX_VALUE;
2149       } else {
2150         comp = new MinFloatComp();
2151         this.nullCompVal = Float.MAX_VALUE;
2152       }
2153 
2154       if(needsScores) {
2155         this.scores = new float[size];
2156         if(nullPolicy == CollapsingPostFilter.NULL_POLICY_EXPAND) {
2157           nullScores = new FloatArrayList();
2158         }
2159       }
2160     }
2161 
2162     public void setNextReader(LeafReaderContext context) throws IOException {
2163       this.minMaxVals = DocValues.getNumeric(context.reader(), this.field);
2164     }
2165 
2166     public void collapse(int collapseKey, int contextDoc, int globalDoc) throws IOException {
2167 
2168       // Check to see if we have documents boosted by the QueryElevationComponent
2169       if(boosts && mergeBoost.boost(globalDoc)) {
2170         boostDocs.add(globalDoc);
2171         boostKeys.add(collapseKey);
2172         return;
2173       }
2174 
2175       int minMaxVal = (int) minMaxVals.get(contextDoc);
2176       float currentVal = Float.intBitsToFloat(minMaxVal);
2177 
2178       if(collapseKey != nullValue) {
2179         final int idx;
2180         if((idx = cmap.indexOf(collapseKey)) >= 0) {
2181           int pointer = cmap.indexGet(idx);
2182           if(comp.test(currentVal, testValues[pointer])) {
2183             testValues[pointer] = currentVal;
2184             docs[pointer] = globalDoc;
2185             if(needsScores) {
2186               scores[pointer] = scorer.score();
2187             }
2188           }
2189         } else {
2190           ++index;
2191           cmap.put(collapseKey, index);
2192           if(index == testValues.length) {
2193             testValues = ArrayUtil.grow(testValues);
2194             docs = ArrayUtil.grow(docs);
2195             if(needsScores) {
2196               scores = ArrayUtil.grow(scores);
2197             }
2198           }
2199 
2200           testValues[index] = currentVal;
2201           docs[index] = globalDoc;
2202           if(needsScores) {
2203             scores[index] = scorer.score();
2204           }
2205         }
2206       } else if(this.nullPolicy == CollapsingPostFilter.NULL_POLICY_COLLAPSE) {
2207         if(comp.test(currentVal, nullCompVal)) {
2208           nullCompVal = currentVal;
2209           nullDoc = globalDoc;
2210           if(needsScores) {
2211             nullScore = scorer.score();
2212           }
2213         }
2214       } else if(this.nullPolicy == CollapsingPostFilter.NULL_POLICY_EXPAND) {
2215         this.collapsedSet.set(globalDoc);
2216         if(needsScores) {
2217           nullScores.add(scorer.score());
2218         }
2219       }
2220     }
2221   }
2222 
2223   /*
2224    *  Strategy for collapsing on a 32 bit numeric field and selecting the group head based
2225    *  on the min/max value of a Value Source Function.
2226    */
2227   private static class IntValueSourceStrategy extends IntFieldValueStrategy {
2228 
2229     private FloatCompare comp;
2230     private float[] testValues;
2231     private float nullCompVal;
2232 
2233     private ValueSource valueSource;
2234     private FunctionValues functionValues;
2235     private Map rcontext;
2236     private final CollapseScore collapseScore = new CollapseScore();
2237     private final boolean cscore;
2238     private float score;
2239     private int index=-1;
2240 
2241     public IntValueSourceStrategy(int maxDoc,
2242                                   int size,
2243                                   String collapseField,
2244                                   int nullValue,
2245                                   int nullPolicy,
2246                                   GroupHeadSelector groupHeadSelector,
2247                                   boolean needsScores,
2248                                   IntIntHashMap boostDocs,
2249                                   FunctionQuery funcQuery,
2250                                   IndexSearcher searcher) throws IOException {
2251 
2252       super(maxDoc, size, collapseField, nullValue, nullPolicy, needsScores, boostDocs);
2253 
2254       this.testValues = new float[size];
2255 
2256       this.valueSource = funcQuery.getValueSource();
2257       this.rcontext = ValueSource.newContext(searcher);
2258 
2259       assert GroupHeadSelectorType.MIN_MAX.contains(groupHeadSelector.type);
2260       
2261       if (GroupHeadSelectorType.MAX.equals(groupHeadSelector.type)) {
2262         this.nullCompVal = -Float.MAX_VALUE;
2263         comp = new MaxFloatComp();
2264       } else {
2265         this.nullCompVal = Float.MAX_VALUE;
2266         comp = new MinFloatComp();
2267       }
2268 
2269       this.cscore = collapseScore.setupIfNeeded(groupHeadSelector, rcontext);
2270 
2271       if(needsScores) {
2272         this.scores = new float[size];
2273         if(nullPolicy == CollapsingPostFilter.NULL_POLICY_EXPAND) {
2274           nullScores = new FloatArrayList();
2275         }
2276       }
2277     }
2278 
2279     public void setNextReader(LeafReaderContext context) throws IOException {
2280       functionValues = this.valueSource.getValues(rcontext, context);
2281     }
2282 
2283     public void collapse(int collapseKey, int contextDoc, int globalDoc) throws IOException {
2284 
2285       // Check to see if we have documents boosted by the QueryElevationComponent
2286       if(boosts && mergeBoost.boost(globalDoc)) {
2287         boostDocs.add(globalDoc);
2288         boostKeys.add(collapseKey);
2289         return;
2290       }
2291 
2292       if(needsScores || cscore) {
2293         this.score = scorer.score();
2294         this.collapseScore.score = score;
2295       }
2296 
2297       float currentVal = functionValues.floatVal(contextDoc);
2298 
2299       if(collapseKey != nullValue) {
2300         final int idx;
2301         if((idx = cmap.indexOf(collapseKey)) >= 0) {
2302           int pointer = cmap.indexGet(idx);
2303           if(comp.test(currentVal, testValues[pointer])) {
2304             testValues[pointer] = currentVal;
2305             docs[pointer] = globalDoc;
2306             if(needsScores){
2307               scores[pointer] = score;
2308             }
2309           }
2310         } else {
2311           ++index;
2312           cmap.put(collapseKey, index);
2313           if(index == testValues.length) {
2314             testValues = ArrayUtil.grow(testValues);
2315             docs = ArrayUtil.grow(docs);
2316             if(needsScores) {
2317               scores = ArrayUtil.grow(scores);
2318             }
2319           }
2320           docs[index] = globalDoc;
2321           testValues[index] = currentVal;
2322           if(needsScores) {
2323             scores[index] = score;
2324           }
2325         }
2326       } else if(this.nullPolicy == CollapsingPostFilter.NULL_POLICY_COLLAPSE) {
2327         if(comp.test(currentVal, nullCompVal)) {
2328           nullCompVal = currentVal;
2329           nullDoc = globalDoc;
2330           if(needsScores) {
2331             nullScore = scorer.score();
2332           }
2333         }
2334       } else if(this.nullPolicy == CollapsingPostFilter.NULL_POLICY_EXPAND) {
2335         this.collapsedSet.set(globalDoc);
2336         if(needsScores) {
2337           nullScores.add(scorer.score());
2338         }
2339       }
2340     }
2341   }
2342 
2343   
2344   /*
2345    * Strategy for collapsing on a 32 bit numeric field and using the first document according 
2346    * to a complex sort as the group head
2347    */
2348   private static class IntSortSpecStrategy extends IntFieldValueStrategy {
2349 
2350     private final SortFieldsCompare compareState;
2351     private final SortSpec sortSpec;
2352     private final Sort sort;
2353     
2354     private float score;
2355     private int index=-1;
2356 
2357     public IntSortSpecStrategy(int maxDoc,
2358                                int size,
2359                                String collapseField,
2360                                int nullValue,
2361                                int nullPolicy,
2362                                GroupHeadSelector groupHeadSelector,
2363                                boolean needsScores,
2364                                IntIntHashMap boostDocs,
2365                                SortSpec sortSpec,
2366                                IndexSearcher searcher) throws IOException {
2367       
2368       super(maxDoc, size, collapseField, nullValue, nullPolicy, needsScores, boostDocs);
2369 
2370       assert GroupHeadSelectorType.SORT.equals(groupHeadSelector.type);
2371 
2372       this.sortSpec = sortSpec;
2373       this.sort = rewriteSort(sortSpec, searcher);
2374       this.compareState = new SortFieldsCompare(sort.getSort(), size);
2375 
2376       if(needsScores) {
2377         this.scores = new float[size];
2378         if(nullPolicy == CollapsingPostFilter.NULL_POLICY_EXPAND) {
2379           nullScores = new FloatArrayList();
2380         }
2381       }
2382     }
2383 
2384     @Override
2385     public void setNextReader(LeafReaderContext context) throws IOException {
2386       compareState.setNextReader(context);
2387     }
2388 
2389     @Override
2390     public void setScorer(Scorer s) {
2391       super.setScorer(s);
2392       this.compareState.setScorer(s);
2393     }
2394 
2395     public void collapse(int collapseKey, int contextDoc, int globalDoc) throws IOException {
2396 
2397       // Check to see if we have documents boosted by the QueryElevationComponent
2398       if(boosts && mergeBoost.boost(globalDoc)) {
2399         boostDocs.add(globalDoc);
2400         boostKeys.add(collapseKey);
2401         return;
2402       }
2403 
2404       if (needsScores) {
2405         this.score = scorer.score();
2406       }
2407 
2408       if (collapseKey != nullValue) {
2409         final int idx;
2410         if ((idx = cmap.indexOf(collapseKey)) >= 0) {
2411           // we've seen this collapseKey before, test to see if it's a new group leader
2412           int pointer = cmap.indexGet(idx);
2413           if (compareState.testAndSetGroupValues(pointer, contextDoc)) {
2414             docs[pointer] = globalDoc;
2415             if (needsScores) {
2416               scores[pointer] = score;
2417             }
2418           }
2419         } else {
2420           // we've never seen this collapseKey before, treat it as group head for now
2421           ++index;
2422           cmap.put(collapseKey, index);
2423           if (index == docs.length) {
2424             docs = ArrayUtil.grow(docs);
2425             compareState.grow(docs.length);
2426             if(needsScores) {
2427               scores = ArrayUtil.grow(scores);
2428             }
2429           }
2430           docs[index] = globalDoc;
2431           compareState.setGroupValues(index, contextDoc);
2432           if(needsScores) {
2433             scores[index] = score;
2434           }
2435         }
2436       } else if(this.nullPolicy == CollapsingPostFilter.NULL_POLICY_COLLAPSE) {
2437         if (-1 == nullDoc) {
2438           // we've never seen a doc with null collapse key yet, treat it as the null group head for now
2439           compareState.setNullGroupValues(contextDoc);
2440           nullDoc = globalDoc;
2441           if (needsScores) {
2442             nullScore = score;
2443           }
2444         } else {
2445           // test this doc to see if it's the new null leader
2446           if (compareState.testAndSetNullGroupValues(contextDoc)) {
2447             nullDoc = globalDoc;
2448             if (needsScores) {
2449               nullScore = score;
2450             }
2451           }
2452         }
2453       } else if(this.nullPolicy == CollapsingPostFilter.NULL_POLICY_EXPAND) {
2454         this.collapsedSet.set(globalDoc);
2455         if (needsScores) {
2456           nullScores.add(score);
2457         }
2458       }
2459     }
2460   }
2461 
2462   static class MergeBoost {
2463 
2464     private int[] boostDocs;
2465     private int index = 0;
2466 
2467     public MergeBoost(int[] boostDocs) {
2468       this.boostDocs = boostDocs;
2469     }
2470 
2471     public void reset() {
2472       this.index = 0;
2473     }
2474 
2475     public boolean boost(int globalDoc) {
2476       if(index == Integer.MIN_VALUE) {
2477         return false;
2478       } else {
2479         while(true) {
2480           if(index >= boostDocs.length) {
2481             index = Integer.MIN_VALUE;
2482             return false;
2483           } else {
2484             int comp = boostDocs[index];
2485             if(comp == globalDoc) {
2486               ++index;
2487               return true;
2488             } else if(comp < globalDoc) {
2489               ++index;
2490             } else {
2491               return false;
2492             }
2493           }
2494         }
2495       }
2496     }
2497   }
2498 
2499   /** 
2500    * Class for comparing documents according to a list of SortField clauses and 
2501    * tracking the groupHeadLeaders and their sort values.  groups will be identified 
2502    * by int "contextKey values, which may either be (encoded) 32bit numeric values, or 
2503    * ordinal values for Strings -- this class doesn't care, and doesn't assume any special 
2504    * meaning.
2505    */
2506   private static class SortFieldsCompare {
2507     final private int numClauses;
2508     final private SortField[] sorts;
2509     final private int[] reverseMul;
2510     final private FieldComparator[] fieldComparators;
2511     final private LeafFieldComparator[] leafFieldComparators;
2512 
2513     private Object[][] groupHeadValues; // growable
2514     final private Object[] nullGroupValues;
2515     
2516     /**
2517      * Constructs an instance based on the the (raw, un-rewritten) SortFields to be used, 
2518      * and an initial number of expected groups (will grow as needed).
2519      */
2520     public SortFieldsCompare(SortField[] sorts, int initNumGroups) throws IOException {
2521       this.sorts = sorts;
2522       numClauses = sorts.length;
2523       fieldComparators = new FieldComparator[numClauses];
2524       leafFieldComparators = new LeafFieldComparator[numClauses];
2525       reverseMul = new int[numClauses];
2526       for (int clause = 0; clause < numClauses; clause++) {
2527         SortField sf = sorts[clause];
2528         // we only need one slot for every comparator
2529         fieldComparators[clause] = sf.getComparator(1, clause);
2530         reverseMul[clause] = sf.getReverse() ? -1 : 1;
2531       }
2532       groupHeadValues = new Object[initNumGroups][];
2533       nullGroupValues = new Object[numClauses];
2534     }
2535 
2536     public void setNextReader(LeafReaderContext context) throws IOException {
2537       for (int clause = 0; clause < numClauses; clause++) {
2538         leafFieldComparators[clause] = fieldComparators[clause].getLeafComparator(context);
2539       }
2540     }
2541     public void setScorer(Scorer s) {
2542       for (int clause = 0; clause < numClauses; clause++) {
2543         leafFieldComparators[clause].setScorer(s);
2544       }
2545     }
2546     
2547     // LUCENE-6808 workarround
2548     private static Object cloneIfBytesRef(Object val) {
2549       if (val instanceof BytesRef) {
2550         return BytesRef.deepCopyOf((BytesRef) val);
2551       }
2552       return val;
2553     }
2554 
2555     /**
2556      * Returns the current SortField values for the specified collapseKey.  
2557      * If this collapseKey has never been seen before, then an array of null values is inited 
2558      * and tracked so that the caller may update it if needed.
2559      */
2560     private Object[] getOrInitGroupHeadValues(int collapseKey) {
2561       Object[] values = groupHeadValues[collapseKey];
2562       if (null == values) {
2563         values = new Object[numClauses];
2564         groupHeadValues[collapseKey] = values;
2565       }
2566       return values;
2567     }
2568       
2569     /**
2570      * Records the SortField values for the specified contextDoc as the "best" values 
2571      * for the group identified by the specified collapseKey. 
2572      *
2573      * Should be called the first time a contextKey is encountered.
2574      */
2575     public void setGroupValues(int collapseKey, int contextDoc) throws IOException {
2576       assert 0 <= collapseKey : "negative collapseKey";
2577       assert collapseKey < groupHeadValues.length : "collapseKey too big -- need to grow array?";
2578       setGroupValues(getOrInitGroupHeadValues(collapseKey), contextDoc);
2579     }
2580     
2581     /**
2582      * Records the SortField values for the specified contextDoc as the "best" values 
2583      * for the null group.
2584      *
2585      * Should be calledthe first time a doc in the null group is encountered
2586      */
2587     public void setNullGroupValues(int contextDoc) throws IOException {
2588       setGroupValues(nullGroupValues, contextDoc);
2589     }
2590     
2591     /**
2592      * Records the SortField values for the specified contextDoc into the 
2593      * values array provided by the caller.
2594      */
2595     private void setGroupValues(Object[] values, int contextDoc) throws IOException {
2596       for (int clause = 0; clause < numClauses; clause++) {
2597         leafFieldComparators[clause].copy(0, contextDoc);
2598         values[clause] = cloneIfBytesRef(fieldComparators[clause].value(0));
2599       }
2600     }
2601 
2602     /**
2603      * Compares the SortField values of the specified contextDoc with the existing group head 
2604      * values for the group identified by the specified collapseKey, and overwrites them
2605      * (and returns true) if this document should become the new group head in accordance 
2606      * with the SortFields
2607      * (otherwise returns false)
2608      */
2609     public boolean testAndSetGroupValues(int collapseKey, int contextDoc) throws IOException {
2610       assert 0 <= collapseKey : "negative collapseKey";
2611       assert collapseKey < groupHeadValues.length : "collapseKey too big -- need to grow array?";
2612       return testAndSetGroupValues(getOrInitGroupHeadValues(collapseKey), contextDoc);
2613     }
2614     
2615     /**
2616      * Compares the SortField values of the specified contextDoc with the existing group head 
2617      * values for the null group, and overwrites them (and returns true) if this document 
2618      * should become the new group head in accordance with the SortFields. 
2619      * (otherwise returns false)
2620      */
2621     public boolean testAndSetNullGroupValues(int contextDoc) throws IOException {
2622       return testAndSetGroupValues(nullGroupValues, contextDoc);
2623     }
2624 
2625     /**
2626      * Compares the SortField values of the specified contextDoc with the existing values
2627      * array, and overwrites them (and returns true) if this document is the new group head in 
2628      * accordance with the SortFields.
2629      * (otherwise returns false)
2630      */
2631     private boolean testAndSetGroupValues(Object[] values, int contextDoc) throws IOException {
2632       Object[] stash = new Object[numClauses];
2633       int lastCompare = 0;
2634       int testClause = 0;
2635       for (/* testClause */; testClause < numClauses; testClause++) {
2636         leafFieldComparators[testClause].copy(0, contextDoc);
2637         FieldComparator fcomp = fieldComparators[testClause];
2638         stash[testClause] = cloneIfBytesRef(fcomp.value(0));
2639         lastCompare = reverseMul[testClause] * fcomp.compareValues(stash[testClause], values[testClause]);
2640         
2641         if (0 != lastCompare) {
2642           // no need to keep checking additional clauses
2643           break;
2644         }
2645       }
2646 
2647       if (0 <= lastCompare) {
2648         // we're either not competitive, or we're completley tied with another doc that's already group head
2649         // that's already been selected
2650         return false;
2651       } // else...
2652       
2653       // this doc is our new group head, we've already read some of the values into our stash
2654       testClause++;
2655       System.arraycopy(stash, 0, values, 0, testClause);
2656       // read the remaining values we didn't need to test
2657       for (int copyClause = testClause; copyClause < numClauses; copyClause++) {
2658         leafFieldComparators[copyClause].copy(0, contextDoc);
2659         values[copyClause] = cloneIfBytesRef(fieldComparators[copyClause].value(0));
2660       }
2661       return true;
2662     }
2663 
2664     /**
2665      * Grows all internal arrays to the specified minSize
2666      */
2667     public void grow(int minSize) {
2668       groupHeadValues = ArrayUtil.grow(groupHeadValues, minSize);
2669     }
2670   }
2671     
2672   private static interface IntCompare {
2673     public boolean test(int i1, int i2);
2674   }
2675 
2676   private static interface FloatCompare {
2677     public boolean test(float i1, float i2);
2678   }
2679 
2680   private static interface LongCompare {
2681     public boolean test(long i1, long i2);
2682   }
2683 
2684   private static class MaxIntComp implements IntCompare {
2685     public boolean test(int i1, int i2) {
2686       return i1 > i2;
2687     }
2688   }
2689 
2690   private static class MinIntComp implements IntCompare {
2691     public boolean test(int i1, int i2) {
2692       return i1 < i2;
2693     }
2694   }
2695 
2696   private static class MaxFloatComp implements FloatCompare {
2697     public boolean test(float i1, float i2) {
2698       return i1 > i2;
2699     }
2700   }
2701 
2702   private static class MinFloatComp implements FloatCompare {
2703     public boolean test(float i1, float i2) {
2704       return i1 < i2;
2705     }
2706   }
2707 
2708   private static class MaxLongComp implements LongCompare {
2709     public boolean test(long i1, long i2) {
2710       return i1 > i2;
2711     }
2712   }
2713 
2714   private static class MinLongComp implements LongCompare {
2715     public boolean test(long i1, long i2) {
2716       return i1 < i2;
2717     }
2718   }
2719 
2720   /** returns the number of arguments that are non null */
2721   private static final int numNotNull(final Object... args) {
2722     int r = 0;
2723     for (final Object o : args) {
2724       if (null != o) {
2725         r++;
2726       }
2727     }
2728     return r;
2729   }
2730 
2731   /**
2732    * Helper method for rewriting the Sort associated with a SortSpec.  
2733    * Handles the special case default of relevancy sort (ie: a SortSpec w/null Sort object)
2734    */
2735   public static Sort rewriteSort(SortSpec sortSpec, IndexSearcher searcher) throws IOException {
2736     assert null != sortSpec : "SortSpec must not be null";
2737     assert null != searcher : "Searcher must not be null";
2738     Sort orig = sortSpec.getSort();
2739     if (null == orig) {
2740       orig = Sort.RELEVANCE;
2741     }
2742     return orig.rewrite(searcher);
2743   }
2744 }